a977c3
@@ -36,9 +36,10 @@
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.ql.stats.StatsUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -55,7 +56,6 @@
 import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
-import org.apache.hadoop.hive.ql.exec.UDTFOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.io.ContentSummaryInputFormat;
@@ -171,7 +171,7 @@
private boolean checkThreshold(FetchData data, int limit, ParseContext pctx) thr
         return true;
       }
     }
-    return data.isDataLengthWitInThreshold(pctx, threshold);
+    return data.isDataLengthWithInThreshold(pctx, threshold);
   }
 
   // all we can handle is LimitOperator, FilterOperator SelectOperator and final FS
@@ -321,6 +321,12 @@
private boolean isConvertible(FetchData fetch, Operator<?> operator, Set<Operato
     return true;
   }
 
+  enum Status {
+    PASS,
+    FAIL,
+    UNAVAILABLE
+  }
+
   private class FetchData {
 
     // source table scan
@@ -417,7 +423,7 @@
private ListSinkOperator completed(ParseContext pctx, FetchWork work) {
       return replaceFSwithLS(fileSink, work.getSerializationNullFormat());
     }
 
-    private boolean isDataLengthWitInThreshold(ParseContext pctx, final long threshold)
+    private boolean isDataLengthWithInThreshold(ParseContext pctx, final long threshold)
         throws Exception {
       if (splitSample != null && splitSample.getTotalLength() != null) {
         if (LOG.isDebugEnabled()) {
@@ -426,74 +432,115 @@
private boolean isDataLengthWitInThreshold(ParseContext pctx, final long thresho
         return (threshold - splitSample.getTotalLength()) > 0;
       }
 
-      final JobConf jobConf = new JobConf(pctx.getConf());
-      Utilities.setColumnNameList(jobConf, scanOp, true);
-      Utilities.setColumnTypeList(jobConf, scanOp, true);
-      HiveStorageHandler handler = table.getStorageHandler();
-      if (handler instanceof InputEstimator) {
-        InputEstimator estimator = (InputEstimator) handler;
-        TableDesc tableDesc = Utilities.getTableDesc(table);
-        PlanUtils.configureInputJobPropertiesForStorageHandler(tableDesc);
-        Utilities.copyTableJobPropertiesToConf(tableDesc, jobConf);
-        long len = estimator.estimate(jobConf, scanOp, threshold).getTotalLength();
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Threshold " + len + " exceeded for pseudoMR mode");
+      Status status = checkThresholdWithMetastoreStats(table, partsList, threshold);
+      if (status.equals(Status.PASS)) {
+        return true;
+      } else if (status.equals(Status.FAIL)) {
+        return false;
+      } else {
+        LOG.info("Cannot fetch stats from metastore for table: {}. Falling back to filesystem scan..",
+          table.getCompleteName());
+        // metastore stats is unavailable, fallback to old way
+        final JobConf jobConf = new JobConf(pctx.getConf());
+        Utilities.setColumnNameList(jobConf, scanOp, true);
+        Utilities.setColumnTypeList(jobConf, scanOp, true);
+        HiveStorageHandler handler = table.getStorageHandler();
+        if (handler instanceof InputEstimator) {
+          InputEstimator estimator = (InputEstimator) handler;
+          TableDesc tableDesc = Utilities.getTableDesc(table);
+          PlanUtils.configureInputJobPropertiesForStorageHandler(tableDesc);
+          Utilities.copyTableJobPropertiesToConf(tableDesc, jobConf);
+          long len = estimator.estimate(jobConf, scanOp, threshold).getTotalLength();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Threshold " + len + " exceeded for pseudoMR mode");
+          }
+          return (threshold - len) > 0;
         }
-        return (threshold - len) > 0;
-      }
-      if (table.isNonNative()) {
-        return true; // nothing can be done
-      }
-      if (!table.isPartitioned()) {
-        long len = getPathLength(jobConf, table.getPath(), table.getInputFormatClass(), threshold);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Threshold " + len + " exceeded for pseudoMR mode");
+        if (table.isNonNative()) {
+          return true; // nothing can be done
         }
-        return (threshold - len) > 0;
-      }
-      final AtomicLong total = new AtomicLong(0);
-      //TODO: use common thread pool later?
-      int threadCount = HiveConf.getIntVar(pctx.getConf(),
+        if (!table.isPartitioned()) {
+          long len = getPathLength(jobConf, table.getPath(), table.getInputFormatClass(), threshold);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Threshold " + len + " exceeded for pseudoMR mode");
+          }
+          return (threshold - len) > 0;
+        }
+        final AtomicLong total = new AtomicLong(0);
+        //TODO: use common thread pool later?
+        int threadCount = HiveConf.getIntVar(pctx.getConf(),
           HiveConf.ConfVars.HIVE_STATS_GATHER_NUM_THREADS);
-      final ExecutorService pool = (threadCount > 0) ?
+        final ExecutorService pool = (threadCount > 0) ?
           Executors.newFixedThreadPool(threadCount,
-              new ThreadFactoryBuilder()
-                  .setDaemon(true)
-                  .setNameFormat("SimpleFetchOptimizer-FileLength-%d").build()) : null;
-      try {
-        List<Future> futures = Lists.newLinkedList();
-        for (final Partition partition : partsList.getNotDeniedPartns()) {
-          final Path path = partition.getDataLocation();
+            new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setNameFormat("SimpleFetchOptimizer-FileLength-%d").build()) : null;
+        try {
+          List<Future> futures = Lists.newLinkedList();
+          for (final Partition partition : partsList.getNotDeniedPartns()) {
+            final Path path = partition.getDataLocation();
+            if (pool != null) {
+              futures.add(pool.submit(new Callable<Long>() {
+                @Override
+                public Long call() throws Exception {
+                  long len = getPathLength(jobConf, path, partition.getInputFormatClass(), threshold);
+                  LOG.trace(path + ", length=" + len);
+                  return total.addAndGet(len);
+                }
+              }));
+            } else {
+              total.addAndGet(getPathLength(jobConf, path, partition.getInputFormatClass(), threshold));
+            }
+          }
           if (pool != null) {
-            futures.add(pool.submit(new Callable<Long>() {
-              @Override
-              public Long call() throws Exception {
-                long len = getPathLength(jobConf, path, partition.getInputFormatClass(), threshold);
-                LOG.trace(path  + ", length=" + len);
-                return total.addAndGet(len);
+            pool.shutdown();
+            for (Future<Long> future : futures) {
+              long totalLen = future.get();
+              if ((threshold - totalLen) <= 0) {
+                // early exit, as getting file lengths can be expensive in object stores.
+                return false;
               }
-            }));
-          } else {
-            total.addAndGet(getPathLength(jobConf, path, partition.getInputFormatClass(), threshold));
-          }
-        }
-        if (pool != null) {
-          pool.shutdown();
-          for (Future<Long> future : futures) {
-            long totalLen = future.get();
-            if ((threshold - totalLen) <= 0) {
-              // early exit, as getting file lengths can be expensive in object stores.
-              return false;
             }
           }
+          return (threshold - total.get()) >= 0;
+        } finally {
+          LOG.info("Data set size=" + total.get() + ", threshold=" + threshold);
+          if (pool != null) {
+            pool.shutdownNow();
+          }
         }
-        return (threshold - total.get()) >= 0;
-      } finally {
-        LOG.info("Data set size=" + total.get() + ", threshold=" + threshold);
-        if (pool != null) {
-          pool.shutdownNow();
+      }
+    }
+
+    // This method gets the basic stats from metastore for table/partitions. This will make use of the statistics from
+    // AnnotateWithStatistics optimizer when available. If execution engine is tez or spark, AnnotateWithStatistics
+    // optimization is applied only during physical compilation because of DPP changing the stats. In such case, we
+    // we will get the basic stats from metastore. When statistics is absent in metastore we will use the fallback of
+    // scanning the filesystem to get file lengths.
+    private Status checkThresholdWithMetastoreStats(final Table table, final PrunedPartitionList partsList,
+      final long threshold) {
+      if (table != null && !table.isPartitioned()) {
+        long dataSize = StatsUtils.getTotalSize(table);
+        if (dataSize <= 0) {
+          LOG.warn("Cannot determine basic stats for table: {} from metastore. Falling back.", table.getCompleteName());
+          return Status.UNAVAILABLE;
         }
+
+        return (threshold - dataSize) >= 0 ? Status.PASS : Status.FAIL;
+      } else if (table != null && table.isPartitioned() && partsList != null) {
+        List<Long> dataSizes = StatsUtils.getBasicStatForPartitions(table, partsList.getNotDeniedPartns(),
+          StatsSetupConst.TOTAL_SIZE);
+        long totalDataSize = StatsUtils.getSumIgnoreNegatives(dataSizes);
+        if (totalDataSize <= 0) {
+          LOG.warn("Cannot determine basic stats for partitioned table: {} from metastore. Falling back.",
+            table.getCompleteName());
+          return Status.UNAVAILABLE;
+        }
+
+        return (threshold - totalDataSize) >= 0 ? Status.PASS : Status.FAIL;
       }
+
+      return Status.UNAVAILABLE;
     }
 
     private long getPathLength(JobConf conf, Path path,
